-
Notifications
You must be signed in to change notification settings - Fork 14.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-9366: Change log4j dependency into log4j2 #7898
Conversation
This requires a KIP since the log4j2 config is not compatible with log4j. |
@ijuma No problem. Thank you for your guidance. 😄 Stay tuned! |
Log4J 2.13.0 contains experimental support for some Log4J 1 Configuration flies. See http://logging.apache.org/log4j/2.x/manual/compatibility.html. |
That's awesome, thanks for sharing. |
I just had to deal with configuring filtered logs in Kafka and was shocked to find it uses log4j 1.2.17. log4j v1 has been dead since 2015. 5 years ago it was known that it should not be used for any new designs and applications should migrate to v2, yet still Apache Kafka stuck with that? Wanted to raise a ticket, but there is this PR, which does not seem to be getting merged anywhere... dear lord... |
@akamensky I am sorry to hear that. I am almost done the KIP and will start the discussion next week. If you are interested in this issue, please join in. Have a nice weekend. |
Man, I really wish more Apache projects started this "upgrade" |
@Cricket007 Sorry for the delay. While working on this issue, I found that this upgrade is much more complicated than I first expected; It is related to lots of module dependencies, API changes, test code modification, and providing backward-compatibility for the logging configuration. Anyway, it is almost done. I successfully upgraded the whole project and now working with some race conditions on test suites. I hope I can complete it in a couple of days. 😉 @akamensky |
No worries. I didn't realize the backwards compatible issues either. I was under the impression that slf4j bridges handled that. Personally, I've been using logback successfully for years |
So, I'm actually working on a project that I just started... Can verify (part of) this log4j-slf4j + slf4j-log4j12 bridging definitely works without issue Notice too <dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
</exclusions>
</dependency> |
import org.apache.log4j.{Level, LogManager, Logger} | ||
import org.apache.logging.log4j.{Level, LogManager} | ||
import org.apache.logging.log4j.core.config.Configurator | ||
import org.apache.logging.log4j.core.LoggerContext |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
IMO, why not rewrite against org.slf4j
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Log4jController
provides a dynamic Logger
querying functionality, not logging itself. It is why it uses log4j Logger
s directly.
In contrast, the streams module does not provide those kinds of functionality so it uses slf4j fascade.
gradle/dependencies.gradle
Outdated
@@ -77,7 +77,7 @@ versions += [ | |||
jersey: "2.28", | |||
jmh: "1.21", | |||
hamcrest: "2.1", | |||
log4j: "1.2.17", | |||
log4j: "2.12.1", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Based on the comments, how about 2.13.x
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@Cricket007 Yes, I am now working with 2.13.1
and it seems like good.
a5eadb9
to
5680e9f
Compare
Here is the WIP update. I almost completed the migration into log4j2
I am also tracking down the reason. If you can give me some advice, it will be a great help! 😃 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
First pass
checkstyle/import-control.xml
Outdated
@@ -276,10 +280,7 @@ | |||
<allow pkg="kafka.utils" /> | |||
<allow pkg="org.apache.zookeeper" /> | |||
<allow pkg="org.apache.zookeeper" /> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is Zookeeper duplicated here??
Also, does Zookeeper transitively bring in log4j anywhere?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This duplication has been addressed in this PR.
Since this setting contols only direct imports only, it does not bring log4j
transitively.
import java.util.Objects; | ||
import java.util.TreeMap; | ||
import org.apache.kafka.connect.errors.NotFoundException; | ||
import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems org.apache.kafka.*
imports used to be first
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed. Please have a look at this PR. :)
.map(logger -> new AbstractMap.SimpleEntry<>(logger.getName(), configLevelToMap(logger))) | ||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is the .map()
needed?
.map(logger -> new AbstractMap.SimpleEntry<>(logger.getName(), configLevelToMap(logger))) | |
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); | |
.collect(Collectors.toMap(logger -> logger.getName(), logger -> configLevelToMap(logger)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also, Collectors.toCollection(Treeset::new)
might be useful
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great. I will apply it. :)
throw new NotFoundException("Logger " + namedLogger + " not found."); | ||
Optional<Logger> found = currentLoggers() | ||
.stream() | ||
.filter(logger -> loggerName.equals(logger.getName())) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can logger == null
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No. log4j2 does not allow logger == null
.
import org.apache.kafka.connect.errors.NotFoundException; | ||
import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems these imports used to be first
Objects.requireNonNull(loggerLevels); | ||
|
||
final LoggerContext loggerContext = Configurator.initialize(LoggingResourceTest.class.getName(), | ||
null, new File("src/test/resources/log4j2.properties").getAbsolutePath()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
personally, rather than rely on src/test/resources, I would pull from the classpath... LoggingResourceTest.class.getClassLoader().getResource("log4j2.properties")
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agree. I will have a try.
gradle/dependencies.gradle
Outdated
@@ -78,6 +78,7 @@ versions += [ | |||
jmh: "1.23", | |||
hamcrest: "2.2", | |||
log4j: "1.2.17", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
remove this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not yet. log4j-appender
is still using log4j; log4j2-appender
is under progress. :)
gradle/dependencies.gradle
Outdated
@@ -165,6 +166,10 @@ libs += [ | |||
kafkaStreams_23: "org.apache.kafka:kafka-streams:$versions.kafka_23", | |||
kafkaStreams_24: "org.apache.kafka:kafka-streams:$versions.kafka_24", | |||
log4j: "log4j:log4j:$versions.log4j", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
remove this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ditto 😃
appender.console.type=Console | ||
appender.console.name=STDOUT | ||
appender.console.layout.type=PatternLayout | ||
appender.console.layout.pattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This pattern looks different
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, but the reason is that it follows deleted quickstart/java/src/main/resources/archetype-resources/src/main/resources/log4j.properties
; it has different pattern so I followed it.
import java.util.Locale; | ||
import java.util.Map; | ||
import java.util.Properties; | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: these got rearranged
@dongjinleekr I hadn't realised you were working on this via KAFKA-9366. I was looking at it via KAFKA-1368. You've made more progress than me, so happy for you to take it forward. But there are a couple of issues I noticed in the course of my effort:
|
5680e9f
to
74e09c5
Compare
Here is the update, with rebasing onto the latest trunk. Now all tests run properly. However, it still has a problem: When I ran the test suites individually (e.g.,
To understand these errors, here is a background: To validate the logging messages, Kafka has been used a test Appender (i.e., This approach works well when running individually, but when running in batch, the test suite randomly fails to initialize the I have been working on this issue for the last week could not find any perfect solution. If any of you have some ideas on this, it would be a great help. Thanks in advance. 😃 |
@tombentley Thanks for your valuable comments - absoultely it will be a great help! I am now applying the comments from @Cricket007 so as soon as it is completed I will review your comments again and leave a feedback. 😄 |
I've written test appenders in other projects just to verify contents of log messages, but not really sure the implications of race conditions on that. |
@Cricket007 It seems like we need to consult to log4j mailing list. Okay, I will have a try. |
What is timeline for merging this PR? |
@dongjinleekr Can please submit a KIP for this? We should have a better good idea of the compatibility implications by now, right? |
Oh, I had missed the comment about the errors we are seeing when running the tests. It may be worth upgrading to the latest release in case it has been fixed. |
All // Sorry for being late, I just got out from my last project; I will have a look at this PR this weekend. |
6f4fd96
to
4c24cf2
Compare
Here is the fix. I completed to implement all the features, migrating tests to follow log4j2 API, and rebasing onto the latest trunk, but there is a problem in logging message validation. When I run
However, If I run the test suites And if I run In contrast, if I run the test methods individually, they also work fine: It seems like there is a problem with log4j in forwarding the log message to the appender. (Or is the appender closed before the log message arrives?) But I can't certain; I followed the way log4j2 test suites do, but could not find similar cases in their codebase. I tried to fix this problem for several days but not succeeded. If you have some spare time, could you check out this PR and run the tests on your machine? I am working with Ubuntu 20.04 + OpenJDK 8. I am curious the same tests also fail in the other environments. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Had a 1st pass to the changes above RocksDBTimestampedStoreTest.java
. Will continue to review the remaining code changes. Thanks.
try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KafkaStreams.class); | ||
final KafkaStreams streams = new KafkaStreams(builder.build(), props, supplier, time)) { | ||
try (final LogCaptureContext logCaptureContext = LogCaptureContext.create(); | ||
final KafkaStreams streams = new KafkaStreams(builder.build(), props, supplier, time)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1
streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
Outdated
Show resolved
Hide resolved
streams/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java
Outdated
Show resolved
Hide resolved
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
Show resolved
Hide resolved
streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java
Outdated
Show resolved
Hide resolved
streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java
Outdated
Show resolved
Hide resolved
streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java
Outdated
Show resolved
Hide resolved
.../streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java
Outdated
Show resolved
Hide resolved
rootLogger.appenderRefs = stdout | ||
rootLogger.appenderRef.stdout.ref = STDOUT | ||
|
||
status = error |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@dongjinleekr , I've had a first pass to all the files. Thanks for the work!
In addition to the comments left, some high level comments:
- Please rebase to the latest trunk to fix merge conflict
- I think we should run a system test and Trogdor test to make sure this change won't break existing tests.
Thank you.
try (final LogCaptureContext logCaptureContext = LogCaptureContext.create()) { | ||
logCaptureContext.setLatch(3); | ||
|
||
// re-open store | ||
try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(RocksDBTimestampedStore.class)) { | ||
// prepare store | ||
rocksDBStore.init((StateStoreContext) context, rocksDBStore); | ||
|
||
assertThat(appender.getMessages(), hasItem("Opening store " + DB_NAME + " in regular mode")); | ||
} finally { | ||
rocksDBStore.put(new Bytes("key".getBytes()), "timestamped".getBytes()); | ||
rocksDBStore.close(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why did we remove finally
block for rocksDBStore.close()
? Is there possible that we have resource leak here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The try block with LogCaptureAppender
resource was removed; That's the reason.
try (LogCaptureAppender appender = LogCaptureAppender.createAndRegister(RocksDBTimestampedStore.class)) { | ||
rocksDBStore.init((StateStoreContext) context, rocksDBStore); | ||
|
||
assertThat(appender.getMessages(), hasItem("Opening store " + DB_NAME + " in upgrade mode")); | ||
} finally { | ||
rocksDBStore.close(); | ||
} | ||
rocksDBStore.init((StateStoreContext) context, rocksDBStore); | ||
assertThat(logCaptureContext.getMessages(), hasItem("INFO Opening store " + DB_NAME + " in upgrade mode ")); | ||
rocksDBStore.close(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto: potential resource leak?
Is there any progress on it? According to this: https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.2.0 , KIP should be completed by March 2 and Feature freeze is on March 16. |
40ec27e
to
5d2fae2
Compare
@showuon Here is the update, rebasing onto the latest trunk. 🙇 |
Hi @dongjinleekr would you consider this patch to fix the compilation error
|
5d2fae2
to
cd8f493
Compare
Rebased onto the latest trunk. cc/ @edoardocomar |
@dongjinleekr Thanks for working on this PR. Is there a timeline on when will this be merged? As per this doc https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.2.0 is it pushed to Kafka 3.3? |
1. Update Dependency Configuration (except log4j-appender, shell, and tools) 1. Change log4j into log4j2 (1.2.7 → 2.14.1) 2. Change log4j dependency into log4j-api + log4j-core 3. Change slf4j-log4j12 dependency into log4j-slf4j-impl 4. Add log4j-1.2-api dependency for backward-compatibility 2. Update core 1. Update Log4jController to use log4j2. (PlaintextAdminIntegrationTest also changed for API change.) 2. Test logging config changed: core/src/test/resources/log4j.properties → core/src/test/resources/log4j2.properties 3. Logging test class changed: LogCaptureAppender → LogCaptureContext, LoggingUtil 4. Enable ignored tests in PlaintextAdminIntegrationTest 3. Update client 1. Test logging config changed: client/src/test/resources/log4j.properties → client/src/test/resources/log4j2.properties 3. Update connect:mirror 1. Test logging config changed: connect/mirror/src/test/resources/log4j.properties → connect/mirror/src/test/resources/log4j2.properties 4. Update connect:runtime 1. Update LoggingResource to use log4j2. 2. Test logging config changed: connect/runtime/src/test/resources/log4j.properties → connect/runtime/src/test/resources/log4j2.properties 5. Update streams 1. Logging test class changed: LogCaptureAppender → LogCaptureContext 2. Test logging config changed: streams/src/test/resources/log4j.properties → streams/src/test/resources/log4j2.properties 3. Add Archetype log4j2 configuration: streams/quickstart/java/src/main/resources/archetype-resources/src/main/resources/log4j2.properties 6. Update streams:test-utils 1. Test logging config changed: streams/test-utils/src/test/resources/log4j.properties → streams/test-utils/src/test/resources/log4j2.properties 7. Update log4j-appender 1. Add a resource-closing process to KafkaLog4jAppenderTest methods that use real Kafka Produce object. (It prevents the tests hanging up.) 8. Update raft 1. Add log4j deprecation message: raft/bin/test-kraft-server-start.sh 2. Add log4j2 configuration: raft/config/kraft-log4j2.properties 9. Update tests - Update tests/kafkatest/services/kafka/kafka.py - tests/kafkatest/services/kafka/templates/log4j.properties → log4j2.properties - Update tests/kafkatest/services/connect.py - tests/kafkatest/services/templates/connect_log4j.properties → connect_log4j2.properties - Update tests/kafkatest/services/trogdor/trogdor.py - tests/kafkatest/services/trogdor/templates/log4j2.properties → log4j2.properties - Update tests/kafkatest/services/streams.py - Add tests/kafkatest/services/templates/tools_log4j2.properties - Update tests/kafkatest/tests/streams/streams_relational_smoke_test.py - tests/kafkatest/tests/streams/templates/log4j_template.properties → log4j2_template.properties 10. Add log4j deprecation messages to the launcher scripts (except tools) - bin/connect-distributed.sh - bin/connect-mirror-maker.sh - bin/connect-standalone.sh - bin/kafka-server-start.sh - bin/zookeeper-server-start.sh - bin/windows/connect-distributed.bat - bin/windows/connect-standalone.bat - bin/windows/kafka-server-start.bat - bin/windows/zookeeper-server-start.bat 11. Add log4j2 conf properties for broker, and connect launcher scripts - config/log4j2.properties - config/connect-log4j2.properties 12. Trivial Cleanups - Remove redundant method call: Defaults.CompressionType.toString → Defaults.CompressionType - Fix typo: the curent root logger level → the current root logger level
…ForLog4jLogLevelsDoesNotWorkWithInvalidConfigs; see KIP-817
cd8f493
to
b7d7b91
Compare
Rebased onto the latest trunk. @dhruvp-8 Sorry for being late. For compatibility reasons, the adoption of this PR is postponed to 4.0. in 3.x, the reloadlog4j will be used instead. You can find out the custom build, patch with log4j2 here - I released the AK 3.1.0 based one this week and working on 3.2.0 based one now. |
Hi @dongjinleekr , we ran into the issue to run zookeepr and kafka , when we tried to use reload4j instead of log4j in kafka-2.8.1 Package. please find below for more details and could you please help us on how to resolve this issue............? Issue : Currently we are using Kafka-2.8.1 in which it has log4j vulnerabilities reported .
After pointing to reload4j, its failing to run kafka and zookeeper with below errors. zookeeper.log kafka.log log4j:ERROR Could not instantiate class [org.apache.log4j.rolling.RollingFileAppender]. could you please help us if we made the changes to those 2 gradle files , is it enought to make kafka-2.8.1 to work using reload4j............? |
@Indupa The correct fully qualified class name for |
@dongjinleekr We're now accepting changes for 4.0 in trunk. Do you think you can rebase this PR? If not, let us know so someone can complete this work. Thanks. |
What is the ETA for 4.0? I might be able to look at it mid-November. |
We're actively working on it in #17373. Hopefully that will be complete for 4.0.0. |
Sorry for being late. I have been very busy nowadays and just got some free time by this end of the year. Thanks for taking the issue. I will join the PR and follow up for the review & improvements. Let's close this PR and continue on #17373! 😃 |
This PR changes log4j dependency into log4j2.
log4j migrated into log4j2 after its last release of 1.2.17 (May 2012), which is affected by this problem. So, the only way to fix it is by moving log4j dependency into log4j2.
The problem is: the API for setting log level dynamically is different between log4j and log4j2. So, this PR also updates how
Log4jController
works.This PR also fixes a potential problem in
Log4jController#getLogLevel
- what if the root logger's level is null? It may result inNullPointerException
.Committer Checklist (excluded from commit message)